Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-35157][runtime] Sources with watermark alignment get stuck once some subtasks finish #24757

Merged
merged 5 commits into from
Jun 14, 2024

Conversation

elon-X
Copy link
Contributor

@elon-X elon-X commented May 6, 2024

What is the purpose of the change

Sources with watermark alignment get stuck once some subtasks finish, this PR solves this problem.

Brief change log

while some subtasks have been finished, the SourceOperator send Long.MAX_VALUE to SourceCoordinator, and SourceCoordinator checks whether subtasks have been finished before sending the event.

Verifying this change

This change added tests and can be verified as follows:

  • org.apache.flink.streaming.api.operators.SourceOperatorAlignmentTest::testWatermarkAlignmentWhileSubtaskFinished()
  • org.apache.flink.test.streaming.api.datastream.WatermarkAlignmentITCase::testTaskFinishedWithWatermarkAlignmentExecution()

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): no
  • The public API, i.e., is any changed class annotated with @Public(Evolving): no
  • The serializers: no
  • The runtime per-record code paths (performance sensitive): no
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: no
  • The S3 file system connector: no

Documentation

  • Does this pull request introduce a new feature? no

@flinkbot
Copy link
Collaborator

flinkbot commented May 6, 2024

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run azure re-run the last Azure build

@elon-X
Copy link
Contributor Author

elon-X commented May 7, 2024

hi, @1996fanrui would you mind reviewing this for me when you have a moment? Thank you very much!

@1996fanrui 1996fanrui self-requested a review May 10, 2024 08:10
@1996fanrui 1996fanrui self-assigned this May 10, 2024
Copy link
Member

@1996fanrui 1996fanrui left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @elon-X , I'm wondering if "NoMoreSplitsEvent" would be appropriate to check if a subtask is finished.

The Split can be discovered dynamically. IIRC, kafka source subtask won't be FINISHED even if it doesn't have split for Flink Streaming Job. (This subtask may be assigned split after adding new kafka partition)

I prefer Gyula proposed solution: The solution could be to send out a max watermark event once the sources finish or to exclude them from the source coordinator.

We need to find a proper code place(source is definitely finished) to send MaxTimestamp.

I guess

may be a suitable place.

@elon-X
Copy link
Contributor Author

elon-X commented Jun 5, 2024

@1996fanrui I've made some changes based on your suggestions. Please review the changes when you have a chance and let me know if there are any further improvements needed. Thanks!

Copy link
Member

@1996fanrui 1996fanrui left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @elon-X , thanks for the update!

The production code looks good to me, I left some comments for test code, please take a look in your free time, thanks~

@@ -377,4 +424,9 @@ private void assertLatestWatermarkAlignmentEvent(int subtask, long expectedWater
assertThat(events.get(events.size() - 1))
.isEqualTo(new WatermarkAlignmentEvent(expectedWatermark));
}

private void assertHasNoMoreSplits(int subtask, boolean expected) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

method name is assertHasNoMoreSplits, but it triggers signalNoMoreSplits as well.

I think sourceCoordinator.getContext().signalNoMoreSplits(subtask); should be moved out of this method.

Also, assertHasNoMoreSplits is only called once, and the method only has one line. I'm not sure whether we need extract it as a separate method. Caller can call assertThat(sourceCoordinator.getContext().hasNoMoreSplits(subtask)).isEqualTo(expected); directly.

assertLatestWatermarkAlignmentEvent(subtask1, 1042);

// mock noMoreSplits event
assertHasNoMoreSplits(subtask0, true);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure what testWatermarkAlignmentWhileSubtaskFinished wants to test?

IIUC, the test still works even if we don't call assertHasNoMoreSplits(subtask0, true); (remove this line) here, right?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My intention is to add a test case to simulate the sending of the noMoreSplits event, indicating that the SourceCoordinator is working correctly. If we have WatermarkAlignmentITCase, perhaps we can remove testWatermarkAlignmentWhileSubtaskFinished. What do you think?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

My intention is to add a test case to simulate the sending of the noMoreSplits event

Actually, this test is testing the watermark result when the watermark of one subtask is Long.Max, right?

I think this test can be removed.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updated, thanks for your patience.

Copy link
Member

@1996fanrui 1996fanrui left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @elon-X for the update!

LGTM assuming the CI is green.

@1996fanrui
Copy link
Member

@elon-X The CI fails, could you rebase the master branch first? We can follow the CI after rebaseing.

@1996fanrui 1996fanrui merged commit 93b9bdf into apache:master Jun 14, 2024
elon-X added a commit to elon-X/flink that referenced this pull request Jun 16, 2024
elon-X added a commit to elon-X/flink that referenced this pull request Jun 16, 2024
elon-X added a commit to elon-X/flink that referenced this pull request Jun 16, 2024
1996fanrui pushed a commit that referenced this pull request Jun 18, 2024
1996fanrui pushed a commit that referenced this pull request Jun 18, 2024
1996fanrui pushed a commit that referenced this pull request Jun 18, 2024
superdiaodiao pushed a commit to superdiaodiao/flink that referenced this pull request Jul 4, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
3 participants